Dagstermill
This library provides an integration with papermill to allow you to run Jupyter notebooks with Dagster.
Related Guides:
- dagstermill.define_dagstermill_asset
- beta
This API is currently in beta, and may have breaking changes in minor version releases, with behavior changes in patch releases.
Creates a Dagster asset for a Jupyter notebook.
Parameters:
- name (str) – The name for the asset
- notebook_path (str) – Path to the backing notebook
- key_prefix (Optional[Union[str, Sequence[str]]]) – If provided, the asset’s key is the concatenation of the key_prefix and the asset’s name, which defaults to the name of the decorated function. Each item in key_prefix must be a valid name in dagster (ie only contains letters, numbers, and _) and may not contain python reserved keywords.
- ins (Optional[Mapping[str, AssetIn]]) – A dictionary that maps input names to information about the input.
- deps (Optional[Sequence[Union[AssetsDefinition, SourceAsset, AssetKey, str]]]) – The assets that are upstream dependencies, but do not pass an input value to the notebook.
- config_schema (Optional[ConfigSchema) – The configuration schema for the asset’s underlying op. If set, Dagster will check that config provided for the op matches this schema and fail if it does not. If not set, Dagster will accept any config provided for the op.
- metadata (Optional[Dict[str, Any]]) – A dict of metadata entries for the asset.
- required_resource_keys (Optional[Set[str]]) – Set of resource handles required by the notebook.
- description (Optional[str]) – Description of the asset to display in the Dagster UI.
- partitions_def (Optional[PartitionsDefinition]) – Defines the set of partition keys that compose the asset.
- op_tags (Optional[Dict[str, Any]]) – A dictionary of tags for the op that computes the asset. Frameworks may expect and require certain metadata to be attached to a op. Values that are not strings will be json encoded and must meet the criteria that json.loads(json.dumps(value)) == value.
- group_name (Optional[str]) – A string name used to organize multiple assets into groups. If not provided, the name “default” is used.
- resource_defs (Optional[Mapping[str, ResourceDefinition]]) – beta (This parameter is currently in beta, and may have breaking changes in minor version releases, with behavior changes in patch releases.) (Beta) A mapping of resource keys to resource definitions. These resources will be initialized during execution, and can be accessed from the context within the notebook.
- io_manager_key (Optional[str]) – A string key for the IO manager used to store the output notebook. If not provided, the default key output_notebook_io_manager will be used.
- retry_policy (Optional[RetryPolicy]) – The retry policy for the op that computes the asset.
- save_notebook_on_failure (bool) – If True and the notebook fails during execution, the failed notebook will be written to the Dagster storage directory. The location of the file will be printed in the Dagster logs. Defaults to False.
- asset_tags (Optional[Dict[str, Any]]) – A dictionary of tags to apply to the asset.
- non_argument_deps (Optional[Union[Set[AssetKey], Set[str]]]) – Deprecated, use deps instead. Set of asset keys that are upstream dependencies, but do not pass an input to the asset.
Examples:
from dagstermill import define_dagstermill_asset
from dagster import asset, AssetIn, AssetKey
from sklearn import datasets
import pandas as pd
import numpy as np
@asset
def iris_dataset():
sk_iris = datasets.load_iris()
return pd.DataFrame(
data=np.c_[sk_iris["data"], sk_iris["target"]],
columns=sk_iris["feature_names"] + ["target"],
)
iris_kmeans_notebook = define_dagstermill_asset(
name="iris_kmeans_notebook",
notebook_path="/path/to/iris_kmeans.ipynb",
ins={
"iris": AssetIn(key=AssetKey("iris_dataset"))
}
)
- dagstermill.define_dagstermill_op
- beta
This API is currently in beta, and may have breaking changes in minor version releases, with behavior changes in patch releases.
Wrap a Jupyter notebook in a op.
Parameters:
- name (str) – The name of the op.
- notebook_path (str) – Path to the backing notebook.
- ins (Optional[Mapping[str, In]]) – The op’s inputs.
- outs (Optional[Mapping[str, Out]]) – The op’s outputs. Your notebook should call
yield_result()
to yield each of these outputs. - required_resource_keys (Optional[Set[str]]) – The string names of any required resources.
- output_notebook_name – (Optional[str]): If set, will be used as the name of an injected output of type of
BufferedIOBase
that is the file object of the executed notebook (in addition to theAssetMaterialization
that is always created). It allows the downstream ops to access the executed notebook via a file object. - asset_key_prefix (Optional[Union[List[str], str]]) – If set, will be used to prefix the asset keys for materialized notebooks.
- description (Optional[str]) – If set, description used for op.
- tags (Optional[Dict[str, str]]) – If set, additional tags used to annotate op. Dagster uses the tag keys notebook_path and kind, which cannot be overwritten by the user.
- io_manager_key (Optional[str]) – If using output_notebook_name, you can additionally provide a string key for the IO manager used to store the output notebook. If not provided, the default key output_notebook_io_manager will be used.
- save_notebook_on_failure (bool) – If True and the notebook fails during execution, the failed notebook will be written to the Dagster storage directory. The location of the file will be printed in the Dagster logs. Defaults to False.
Returns:
OpDefinition
class
dagstermill.ConfigurableLocalOutputNotebookIOManager- beta
This API is currently in beta, and may have breaking changes in minor version releases, with behavior changes in patch releases.
Built-in IO Manager for handling output notebook.
- dagstermill.get_context
Get a dagstermill execution context for interactive exploration and development.
Parameters:
- op_config (Optional[Any]) – If specified, this value will be made available on the context as its
op_config
property. - resource_defs (Optional[Mapping[str, ResourceDefinition]]) – Specifies resources to provide to context.
- logger_defs (Optional[Mapping[str, LoggerDefinition]]) – Specifies loggers to provide to context.
- run_config (Optional[dict]) – The config dict with which to construct the context.
Returns:
DagstermillExecutionContext
- op_config (Optional[Any]) – If specified, this value will be made available on the context as its
- dagstermill.yield_event
Yield a dagster event directly from notebook code.
When called interactively or in development, returns its input.
Parameters: dagster_event (Union[
dagster.AssetMaterialization
,dagster.ExpectationResult
,dagster.TypeCheck
,dagster.Failure
,dagster.RetryRequested
]) – An event to yield back to Dagster.
- dagstermill.yield_result
Yield a result directly from notebook code.
When called interactively or in development, returns its input.
Parameters:
- value (Any) – The value to yield.
- output_name (Optional[str]) – The name of the result to yield (default:
'result'
).
class
dagstermill.DagstermillExecutionContext- beta
This API is currently in beta, and may have breaking changes in minor version releases, with behavior changes in patch releases.
Dagstermill-specific execution context.
Do not initialize directly: use
dagstermill.get_context()
.property
job_defThe job definition for the context.
This will be a dagstermill-specific shim.
Type:
dagster.JobDefinition
property
job_nameThe name of the executing job.
Type: str
property
logging_tagsThe logging tags for the context.
Type: dict
property
op_configA dynamically-created type whose properties allow access to op-specific config.
Type: collections.namedtuple
property
op_defThe op definition for the context.
In interactive contexts, this may be a dagstermill-specific shim, depending whether an op definition was passed to
dagstermill.get_context
.Type:
dagster.OpDefinition
property
runThe job run for the context.
Type:
dagster.DagsterRun
property
run_configThe run_config for the context.
Type: dict
property
run_idThe run_id for the context.
Type: str
class
dagstermill.DagstermillErrorBase class for errors raised by dagstermill.